Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

opt,sql: use paired-joins with non-covering indexes for left joins #58261

Merged
merged 1 commit into from
Feb 2, 2022

Conversation

sumeerbhola
Copy link
Collaborator

This is done when the left outer/semi/anti join can use a
lookup join. Prior to this, when the non-covering index
could not fully evaluate the filter for left join we could
not generate a lookup join.

With this change:

  • Left outer join becomes a pair of two left outer joins.
  • Left semi join is a pair of inner join followed by left
    semi join.
  • Left anti join is a pair of left outer join followed by
    left anti join.

Informs #55452

Release note (performance improvement): The optimizer can now
generate lookup joins in certain cases for non-covering
indexes, when performing a left outer/semi/anti join.

@sumeerbhola sumeerbhola requested a review from a team as a code owner December 23, 2020 22:14
@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've probably missed some places to change, and this may need more testing.
Also, I need guidance on how to fix the ordering issue: the paired-joins require the first join to preserve the input ordering. Currently I've put a hack in distsql_physical_planner.go to set MaintainOrdering to true, but this should really happen earlier in the optimizer when doing transformations.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @rytaft)

Copy link
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice! Can we remove the ConvertSemiToInnerJoin rule now?

The ordering issue is a bit tricky, since I don't think you want to force a sort on the input -- you just want to make sure that rows originating from the same input row don't get separated.

Also, at the moment, we don't even take into account the need to maintain ordering in our cost model for lookup joins. So I'm not sure you can do much better than the hack you've added (although we could potentially move the hack into the execbuilder). Anything we add in the optimizer to enforce this ordering will probably add some unintended overhead during execution.

@RaduBerinde do you think there is a way to do this cleanly in the optimizer?

Reviewed 15 of 15 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @sumeerbhola)


pkg/sql/opt/xform/join_funcs.go, line 361 at r1 (raw file):

In some cases

I'd make clear that this only applies for semi, anti, and some left joins


pkg/sql/opt/xform/join_funcs.go, line 363 at r1 (raw file):

		// In some cases, this lower-upper pair of joins is further specialized
		// into paired-joins where we refer to the lower as first and upper as
		// second. The paired-joins use a continuation column in the first join.

I think this could use more detail. It would be great if you can update the function comment to describe this case. Might want to separate it into "case 3".


pkg/sql/opt/xform/join_funcs.go, line 455 at r1 (raw file):

			indexJoin.IsSecondJoinInPairedJoiner = true
		}

Is it worth putting the rest of this into a helper function? Seems like this is basically identical to the code in GenerateInvertedJoins.


pkg/sql/opt/xform/join_funcs.go, line 468 at r1 (raw file):

		// lookup join).
		outputCols := indexJoin.Cols.Intersection(indexJoin.Input.Relational().OutputCols)
		if outputCols.SubsetOf(input.Relational().OutputCols) {

[nit] to be consistent with the rest of this function, use inputProps.OutputCols instead of input.Relational().OutputCols.

Copy link
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTR!
Do we have any existing benchmarks with non-covering indices that I can try this on, or some execution benchmarks that I could extend?

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @sumeerbhola)

@rytaft
Copy link
Collaborator

rytaft commented Jan 5, 2021

There are no optimizer-specific benchmarks for this, but I bet you could modify one of the workloads in the workload directory. TPC-DS has a bunch of left joins (although not sure what the status is of that benchmark and if it's runnable). Probably worth asking around in Slack...

Copy link
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @rytaft)


pkg/sql/opt/xform/join_funcs.go, line 361 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…
In some cases

I'd make clear that this only applies for semi, anti, and some left joins

Done


pkg/sql/opt/xform/join_funcs.go, line 363 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

I think this could use more detail. It would be great if you can update the function comment to describe this case. Might want to separate it into "case 3".

Done.


pkg/sql/opt/xform/join_funcs.go, line 455 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

Is it worth putting the rest of this into a helper function? Seems like this is basically identical to the code in GenerateInvertedJoins.

Good point. Done


pkg/sql/opt/xform/join_funcs.go, line 468 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

[nit] to be consistent with the rest of this function, use inputProps.OutputCols instead of input.Relational().OutputCols.

Done.


pkg/sql/opt/xform/join_funcs.go, line 161 at r2 (raw file):

//     and then use a lookup join to retrieve column z. The "index join" (top
//     LookupJoin) will produce columns a,b,c,x,y,z; the lookup columns are just x
//     (the original index join produced a,b,c,x,y).

I changed this comment since it seemed incorrect, though maybe I misunderstood it. PTAL.

Copy link
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 2 of 3 files at r2, 4 of 4 files at r3.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @sumeerbhola)


pkg/sql/opt/xform/join_funcs.go, line 455 at r1 (raw file):

Previously, sumeerbhola wrote…

Good point. Done

Can you call the function from GenerateInvertedJoins too?


pkg/sql/opt/xform/join_funcs.go, line 161 at r2 (raw file):

Previously, sumeerbhola wrote…

I changed this comment since it seemed incorrect, though maybe I misunderstood it. PTAL.

I think this should still say "the lookup columns are just z", but the other changes you made look right to me.

Also might want to change "original index join" to say "original lookup join".


pkg/sql/opt/xform/join_funcs.go, line 174 at r3 (raw file):

//     joins, but paired-joins explicitly know their role in the pair and
//     behave accordingly.
//     For example, using the same tables in the example for case 2:

[nit] add an empty line above this one


pkg/sql/opt/xform/join_funcs.go, line 175 at r3 (raw file):

//     behave accordingly.
//     For example, using the same tables in the example for case 2:
//      SELECT * FROM abc JOIN xyz ON a=y AND b=z

maybe change this to LEFT JOIN so it would actually use paired joins


pkg/sql/opt/xform/join_funcs.go, line 484 at r3 (raw file):

}

// addIndexJoinAsSecondJoin is a helper function used when constructing two

[nit] this should match the name of the function. But I kind of like this name better than the function name... I'd remove the "ToMemo" from the function name, since I think that's implied.

Copy link
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, I need guidance on how to fix the ordering issue: the paired-joins require the first join to preserve the input ordering. Currently I've put a hack in distsql_physical_planner.go to set MaintainOrdering to true, but this should really happen earlier in the optimizer when doing transformations.

The ordering issue is a bit tricky, since I don't think you want to force a sort on the input -- you just want to make sure that rows originating from the same input row don't get separated.

I don't think MaintainOrdering=true will force a sort on the input -- it switches the joinReader to use joinReaderOrderingStrategy that needs to buffer the looked up rows (unlike joinReaderNoOrderingStrategy) so it can produce all the output rows for an input row in-order, which may result in spilling to disk, so can be more expensive.

Also, at the moment, we don't even take into account the need to maintain ordering in our cost model for lookup joins. So I'm not sure you can do much better than the hack you've added (although we could potentially move the hack into the execbuilder). Anything we add in the optimizer to enforce this ordering will probably add some unintended overhead during execution.

@RaduBerinde do you think there is a way to do this cleanly in the optimizer?

This ordering issue is the main open issue. I'd like a cleaner solution.
@RaduBerinde do you have any thoughts on this?

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @rytaft)


pkg/sql/opt/xform/join_funcs.go, line 455 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

Can you call the function from GenerateInvertedJoins too?

It does below. Maybe I misunderstood your comment.


pkg/sql/opt/xform/join_funcs.go, line 161 at r2 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

I think this should still say "the lookup columns are just z", but the other changes you made look right to me.

Also might want to change "original index join" to say "original lookup join".

The term "lookup columns" seems poorly defined. In LookupJoinPrivate there are comments like:
"set of lookup columns retrieved through lookup" (which suggests z), and "lookup columns form a key in the table" (which suggests x), and the JoinReaderSpec comment "(lookup columns is [0])" also suggest x.
I am changing it back to z for now.

And changed to "original lookup join".


pkg/sql/opt/xform/join_funcs.go, line 174 at r3 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

[nit] add an empty line above this one

Done.


pkg/sql/opt/xform/join_funcs.go, line 175 at r3 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

maybe change this to LEFT JOIN so it would actually use paired joins

Done.


pkg/sql/opt/xform/join_funcs.go, line 484 at r3 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

[nit] this should match the name of the function. But I kind of like this name better than the function name... I'd remove the "ToMemo" from the function name, since I think that's implied.

Done. I changed one and forgot to change the other.

Copy link
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think MaintainOrdering=true will force a sort on the input

Right, I agree. What I meant was that I think if you try to maintain ordering from the perspective of the optimizer, it will force a sort on the input. I don't think we have a way for the optimizer to set MaintainOrdering=true unless there is an explicit ordering.

@RaduBerinde might have some insight, but he's out of office until next week.

Reviewed 1 of 1 files at r4.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @sumeerbhola)


pkg/sql/opt/xform/join_funcs.go, line 455 at r1 (raw file):

Previously, sumeerbhola wrote…

It does below. Maybe I misunderstood your comment.

Sorry! Not sure how I missed that.


pkg/sql/opt/xform/join_funcs.go, line 161 at r2 (raw file):

Previously, sumeerbhola wrote…

The term "lookup columns" seems poorly defined. In LookupJoinPrivate there are comments like:
"set of lookup columns retrieved through lookup" (which suggests z), and "lookup columns form a key in the table" (which suggests x), and the JoinReaderSpec comment "(lookup columns is [0])" also suggest x.
I am changing it back to z for now.

And changed to "original lookup join".

Yea, I agree it's confusing. You could also change "lookup columns" to say "the columns retrieved from the primary index". But up to you -- LGTM otherwise.

Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If I understand correctly, the paired joins are guaranteed to be paired 1-to-1 with no data shuffling between the two stages, correct? If that's the case, we can store the continuation column in both the upper and the lower lookup join and have the upper join always require an ordering on the continuation column. The code in lookupOrIndexJoinCanProvideOrdering would be changed to reflect that a lookup join can always provide an ordering on the continuation column. We're lying to distsql that we have an ordering on that column, but that's ok since we don't use ordered synchronizers if there is a single input stream.

I don't think this would be better than what you have here though (seems to introduce more confusion). If we care about any costing differences, we can directly take into account flags like IsFirstJoinInPairedJoiner in the coster.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola)


pkg/sql/opt/ops/relational.opt, line 343 at r4 (raw file):

    # IsFirstJoinInPairedJoiner is true if this is the first join of a
    # paired-joiner used for left joins.
    IsFirstJoinInPairedJoiner bool

[nit] I would use "lower" and "upper" everywhere, since it's not obvious which is first and which is second (depends if you think about the plan tree or the flow of data).

Copy link
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(trying to translate this into something I understand, given my limited understanding of the optimizer code)

If I understand correctly, the paired joins are guaranteed to be paired 1-to-1 with no data shuffling between the two stages, correct?

Yes, we want them to stay paired with no data shuffling. If data is shuffled between the two, the result will be incorrect. The existing if-block in lookupOrIndexJoinCanProvideOrdering

if lookupJoin, ok := expr.(*memo.LookupJoinExpr); ok && lookupJoin.IsSecondJoinInPairedJoiner {
// Can only pass through ordering if the ordering can be provided by the
// child, since we don't want a sort to be interposed between the child
// and this join.
return CanProvide(expr.Child(0).(memo.RelExpr), required)
}
attempts to ensure this, but I'd like to confirm that this is indeed sufficient. There is also the code in CheckExpr for tests.

If that's the case, we can store the continuation column in both the upper and the lower lookup join and have the upper join always require an ordering on the continuation column. The code in lookupOrIndexJoinCanProvideOrdering would be changed to reflect that a lookup join can always provide an ordering on the continuation column.

The continuation column is a bool, and we expect each input to produce a sequence of (false, true*). Actually sorting by it would be incorrect.
If the upper join also provided a continuation column, I am assuming we would then need to add a project to remove it, yes?
Also, I'm unclear on how lookupOrIndexJoinCanProvideOrdering can claim to provide ordering on the continuation column, given that it is just answering true/false to a question it is being asked regarding the OrderingChoice parameter. Who would adjust the OrderingChoice parameter to include the continuation col? And there could be other expressions in the same memo group as the paired-joins that don't provide the continuation column -- I'm not clear how that would work.

We're lying to distsql that we have an ordering on that column, but that's ok since we don't use ordered synchronizers if there is a single input stream.

I am assuming you mean that each stream from a lower joinReader will have a corresponding upper joinReader, so there won't be an OrderedSynchronizer between the two. Correct?
But what about above the upper joinReader, since it is also claiming to provide an ordered continuation column -- couldn't there be an OrderedSynchronizer there to merge streams from multiple upper joinReaders?

I don't think this would be better than what you have here though (seems to introduce more confusion). If we care about any costing differences, we can directly take into account flags like IsFirstJoinInPairedJoiner in the coster.

Are you suggesting that we would change computeIndexLookupJoinCost in coster.go to look at whether memo.LookupJoinExpr has IsFirstJoinInPairedJoiner set, and if so change its costs knowing that it will respect input ordering? I can add a TODO there, for when the optimizer starts taking ordering cost into account for lookup joins. Or if you all already have a cost model in mind, I can add it now.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola)

Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not suggesting that the upper lookup join produces a continuation column. I am suggesting that it knows the ID of the continuation column from the lower join, so that it can always require an ordering on that column from its child (that would happen in lookupOrIndexJoinBuildChildReqOrdering); then the canProvide in the child would know that it can always provide that ordering. I understand that the column isn't sorted in reality, we would just be pretending from the optimizer side. It definitely doesn't fix any core issue (and in fact, we will probably need special code to know that this special ordering can't be enforced with a sort).

In any case, we would still be relying on two things:

We could upgrade the continuation column to an int64 with an increasing "group" index. Then we would actually have a true ordering on this column (and it wouldn't be incorrect to enforce it by sorting). If we mix in a host ID or random value, we could in principle allow for multiple lower joiners to feed in a single upper joiner with an ordered synchronizer. But this feels unnecessary in the current implementation.

I would just go with your current solution in the PR for now. Longer term, we should brainstorm on what the missing concept is. The point of the continuation column is to be able to group the results for a single input row, is this correct? I think we could achieve that in a much more elegant fashion by wrapping the input in an Ordinality and then plumbing through that ordinality column. We would have a true ordering on it and we wouldn't need to create any special columns. Ordinality doesn't get distributed, but if we care about that, we could add a DistributedOrdinality variant that adds a node id or similar to the output value to prevent collisions (we could use that in some other cases as well).

And there could be other expressions in the same memo group as the paired-joins that don't provide the continuation column

Oh.. I assumed that these lookup joins produce the continuation column as an output column (which would mean they can't be in the same group with expressions that don't produce that column). It looks like they don't, which is pretty sketchy. Until they do, we can't do anything like requiring an ordering on that column.

Are you suggesting that we would change computeIndexLookupJoinCost in coster.go to look at whether memo.LookupJoinExpr has IsFirstJoinInPairedJoiner set, and if so change its costs knowing that it will respect input ordering?

Right, you can just add a TODO.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola)


pkg/sql/opt/exec/execbuilder/relational.go, line 1477 at r4 (raw file):

		}
		// Assign the continuation column the next unused value in the map.
		allCols.Set(int(join.ContinuationCol), maxValue+1)

Is it possible for a first paired join to be an anti or semi? Because the column wouldn't be in the map in that case (see the following few lines)

Copy link
Collaborator Author

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTRs!

I'll hold off on merging until we have a solution for #58892

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola)

@rytaft
Copy link
Collaborator

rytaft commented Feb 22, 2021

Should this be merged? #58892 was fixed a while ago. Would be great to get this into the release!

@sumeerbhola
Copy link
Collaborator Author

I need to rework this in light of that fix, but I am unlikely to have time to work on it this week. If you or someone else has time this week, please feel free to take this over.

@rytaft
Copy link
Collaborator

rytaft commented Feb 23, 2021

We discussed during our weekly meeting and it doesn't seem like any of us on the optimizer team will have free cycles this week. If you don't have a chance to get to it, hopefully we can get this in early in the next release cycle.

@tbg tbg added the X-noremind Bots won't notify about PRs with X-noremind label May 6, 2021
@yuzefovich
Copy link
Member

Pinging @cockroachdb/sql-queries in case anyone is curious to push this over the finish line (btw we have fix-it day on Monday :) ).

@sumeerbhola
Copy link
Collaborator Author

@rytaft any possibility of someone picking this up? It mainly needs someone with the latest optimizer knowledge to do a well-thought rebase onto master. I am unlikely to come back to this given my stale knowledge of that code base, and other work.

@rytaft
Copy link
Collaborator

rytaft commented Jan 26, 2022

sorry for the delayed response. I'll take this over and try to push it over the finish line

Copy link
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did the rebase, and all the opt tests are passing now. Let's see if any other tests fail. Otherwise, I think this is ready to go. Does anyone from SQL Queries want to take a quick glance at this to make sure I didn't miss something?

Reviewed 16 of 16 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @sumeerbhola)


pkg/sql/opt/exec/execbuilder/relational.go, line 1477 at r4 (raw file):

Previously, RaduBerinde wrote…

Is it possible for a first paired join to be an anti or semi? Because the column wouldn't be in the map in that case (see the following few lines)

No I think the first join is always either a left or an inner join.


pkg/sql/opt/ops/relational.opt, line 343 at r4 (raw file):

Previously, RaduBerinde wrote…

[nit] I would use "lower" and "upper" everywhere, since it's not obvious which is first and which is second (depends if you think about the plan tree or the flow of data).

I'm going to leave this as-is for now since first / second is already used throughout the codebase and this would be a big change.

@rytaft rytaft requested a review from a team January 28, 2022 23:13
@rytaft rytaft removed the X-noremind Bots won't notify about PRs with X-noremind label Jan 28, 2022
Copy link
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 1 of 1 files at r6.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde and @sumeerbhola)

Copy link
Member

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did a quick pass. Probably it'd be good to get an approval from @RaduBerinde.

Reviewed 15 of 16 files at r5, 1 of 1 files at r6.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde, @rytaft, and @sumeerbhola)


pkg/sql/lookup_join.go, line 69 at r6 (raw file):

	// At most one of is{First,Second}JoinInPairedJoiner can be true.
	// IsFirstJoinInPairedJoiner can be true only if reqOrdering asks the join

Doesn't this comment contradict the TODO added in the distsql_physical_planner?


pkg/sql/opt/exec/execbuilder/relational.go, line 1477 at r4 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

No I think the first join is always either a left or an inner join.

nit: maybe add this observation as a comment.


pkg/sql/opt/xform/join_funcs.go, line 674 at r6 (raw file):

				// The lowerJoinType continues to be LeftJoinOp.
			}
			// We have already set pairedJoins=true for SemiJoin,AntiJoin earlier,

nit: add a space before AntiJoin.


pkg/sql/opt/exec/execbuilder/testdata/lookup_join, line 774 at r6 (raw file):

# Left anti-join with ON filter on non-covering index
query T
EXPLAIN (VERBOSE) SELECT small.c FROM small WHERE NOT EXISTS(SELECT 1 FROM large WHERE small.c = large.b AND large.d < small.a)

nit: this query doesn't seem to match to what is added in the logic tests above.

@rytaft rytaft force-pushed the paired_lookup branch 2 times, most recently from d30ecb9 to 99bd67a Compare January 31, 2022 20:38
Copy link
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTR! Updated.

Reviewed 3 of 3 files at r7, 1 of 1 files at r8.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @RaduBerinde, @sumeerbhola, and @yuzefovich)


pkg/sql/lookup_join.go, line 69 at r6 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

Doesn't this comment contradict the TODO added in the distsql_physical_planner?

Good point. Fixed.


pkg/sql/opt/exec/execbuilder/relational.go, line 1477 at r4 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: maybe add this observation as a comment.

I added an assertion


pkg/sql/opt/xform/join_funcs.go, line 674 at r6 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: add a space before AntiJoin.

Done.


pkg/sql/opt/exec/execbuilder/testdata/lookup_join, line 774 at r6 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: this query doesn't seem to match to what is added in the logic tests above.

Done.

@rytaft rytaft requested a review from a team January 31, 2022 21:50
Copy link
Collaborator

@mgartner mgartner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cool! :lgtm:

Reviewed 11 of 16 files at r5, 1 of 1 files at r6, 3 of 3 files at r7, 1 of 1 files at r8.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @RaduBerinde, @rytaft, @sumeerbhola, and @yuzefovich)


pkg/sql/opt_exec_factory.go, line 698 at r8 (raw file):

	n.columns = pred.cols
	if isFirstJoinInPairedJoiner {
		n.columns = append(n.columns, colinfo.ResultColumn{Name: "cont", Typ: types.Bool})

It seems dangerous to assume the continuation column is always the last column. Is there no way to use the continuation column in the LookupJoinPrivate to make this more robust?


pkg/sql/opt/memo/check_expr.go, line 250 at r8 (raw file):

				}
			default:
				panic(errors.AssertionFailedf("lookup paired-join is paired with %T", t.Input))

nit: you could also add validation for the continuation columm


pkg/sql/opt/ops/relational.opt, line 343 at r4 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

I'm going to leave this as-is for now since first / second is already used throughout the codebase and this would be a big change.

Can you expand these comments to clarify which join is first here which is second?


pkg/sql/opt/ops/relational.opt, line 416 at r8 (raw file):


    # ContinuationCol is the column ID of the continuation column when
    # IsFirstJoinInPairedJoiner is true.

nit: an explanation of this continuation column might be nice here


pkg/sql/opt/xform/join_funcs.go, line 204 at r8 (raw file):

//     where cont is the continuation column used to group together rows that
//     correspond to the same original a,b,c. The second join will evaluate
//     b=z and produce columns a,b,c,x,y,z. A similar approach works for

Might be more clear to say: "The second join will fetch z from the primary index, evaluate b=z, and produce ...". I was confused because I originally thought it meant the second lookup join will have key columns b and z, which wouldn't be possible without an index on z.


pkg/sql/opt/xform/testdata/rules/join, line 2339 at r8 (raw file):

 │    ├── columns: m:1 n:2 a:6 b:7 abcd.rowid:9 continuation:12
 │    ├── key columns: [1] = [6]
 │    ├── fd: (9)-->(6,7,12)

nit: it might be helpful to print first join in paired joiner and the continuation column in the expression tree

This is done when the left outer/semi/anti join can use a
lookup join. Prior to this, when the non-covering index
could not fully evaluate the filter for left join we could
not generate a lookup join.

With this change:
- Left outer join becomes a pair of two left outer joins.
- Left semi join is a pair of inner join followed by left
  semi join.
- Left anti join is a pair of left outer join followed by
  left anti join.

Informs cockroachdb#55452

Release note (performance improvement): The optimizer can now
generate lookup joins in certain cases for non-covering
indexes, when performing a left outer/semi/anti join.
Copy link
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTR!

Reviewed 8 of 8 files at r9.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @mgartner, @sumeerbhola, and @yuzefovich)


pkg/sql/opt_exec_factory.go, line 698 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

It seems dangerous to assume the continuation column is always the last column. Is there no way to use the continuation column in the LookupJoinPrivate to make this more robust?

I don't really see a way to do that here. We already have that logic inside the execbuilder.


pkg/sql/opt/exec/execbuilder/relational.go, line 1477 at r4 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

I added an assertion

Moved the assertion to check_expr


pkg/sql/opt/exec/execbuilder/relational.go, line 1769 at r8 (raw file):

				join.JoinType.String(),
			)
		}

Note that I moved this assertion to check_expr


pkg/sql/opt/memo/check_expr.go, line 250 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

nit: you could also add validation for the continuation columm

Done.


pkg/sql/opt/ops/relational.opt, line 343 at r4 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

Can you expand these comments to clarify which join is first here which is second?

Done.


pkg/sql/opt/ops/relational.opt, line 416 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

nit: an explanation of this continuation column might be nice here

Done.


pkg/sql/opt/xform/join_funcs.go, line 204 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

Might be more clear to say: "The second join will fetch z from the primary index, evaluate b=z, and produce ...". I was confused because I originally thought it meant the second lookup join will have key columns b and z, which wouldn't be possible without an index on z.

Done.


pkg/sql/opt/xform/testdata/rules/join, line 2339 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

nit: it might be helpful to print first join in paired joiner and the continuation column in the expression tree

Done.

Copy link
Collaborator

@rytaft rytaft left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

bors r+

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @mgartner, @sumeerbhola, and @yuzefovich)

@craig craig bot merged commit 88522a3 into cockroachdb:master Feb 2, 2022
@craig
Copy link
Contributor

craig bot commented Feb 2, 2022

Build succeeded:

Copy link
Collaborator

@mgartner mgartner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 8 of 8 files at r9.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale)


pkg/sql/opt/exec/execbuilder/relational.go, line 1769 at r8 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

Note that I moved this assertion to check_expr

I'm not sure it matters, but perviously this assertion would run for release builds, but in check_expr it will only run for test builds.

@rytaft
Copy link
Collaborator

rytaft commented Feb 2, 2022


pkg/sql/opt/exec/execbuilder/relational.go, line 1769 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

I'm not sure it matters, but perviously this assertion would run for release builds, but in check_expr it will only run for test builds.

Yea, I think that there are a lot of things we could assert in the execbuilder that we don't since we rely on them being caught during testing by assertions in check_expr. It seemed more consistent to follow that convention.

Also note that I had just added that assertion as part of this PR, so it was never running before.

@sumeerbhola
Copy link
Collaborator Author

@rytaft Thanks a lot for pushing this over the finish line!

@rytaft
Copy link
Collaborator

rytaft commented Feb 9, 2022

np! Thank you for the PR and for the nudge to finish it!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants